f7add7
@@ -26,6 +26,7 @@
import java.io.OutputStream;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.io.compress.CodecPool;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.CompressionInputStream;
@@ -34,6 +35,7 @@
import org.apache.hadoop.io.compress.Compressor;
 import org.apache.hadoop.io.compress.Decompressor;
 import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.util.ReflectionUtils;
 
 /**
  * Compression related stuff.
@@ -74,24 +76,19 @@
public final class Compression {
   public static enum Algorithm {
     LZO("lzo") {
       // Use base type to avoid compile-time dependencies.
-      private DefaultCodec lzoCodec;
+      private CompressionCodec lzoCodec;
 
       @Override
-      DefaultCodec getCodec() {
+      CompressionCodec getCodec() {
         if (lzoCodec == null) {
           Configuration conf = new Configuration();
           conf.setBoolean("hadoop.native.lib", true);
           try {
             Class externalCodec =
                 ClassLoader.getSystemClassLoader().loadClass("com.hadoop.compression.lzo.LzoCodec");
-            lzoCodec = (DefaultCodec) externalCodec.newInstance();
-            lzoCodec.setConf(conf);
+            lzoCodec = (CompressionCodec) ReflectionUtils.newInstance(externalCodec, conf);
           } catch (ClassNotFoundException e) {
             throw new RuntimeException(e);
-          } catch (InstantiationException e) {
-            throw new RuntimeException(e);
-          } catch (IllegalAccessException e) {
-            throw new RuntimeException(e);
           }
         }
         return lzoCodec;
@@ -156,15 +153,16 @@
public final class Compression {
       this.compressName = name;
     }
 
-    abstract DefaultCodec getCodec();
+    abstract CompressionCodec getCodec();
 
     public InputStream createDecompressionStream(
         InputStream downStream, Decompressor decompressor,
         int downStreamBufferSize) throws IOException {
-      DefaultCodec codec = getCodec();
+      CompressionCodec codec = getCodec();
       // Set the internal buffer size to read from down stream.
       if (downStreamBufferSize > 0) {
-        codec.getConf().setInt("io.file.buffer.size", downStreamBufferSize);
+        Configurable c = (Configurable) codec;
+        c.getConf().setInt("io.file.buffer.size", downStreamBufferSize);
       }
       CompressionInputStream cis =
           codec.createInputStream(downStream, decompressor);
@@ -176,7 +174,7 @@
public final class Compression {
     public OutputStream createCompressionStream(
         OutputStream downStream, Compressor compressor, int downStreamBufferSize)
         throws IOException {
-      DefaultCodec codec = getCodec();
+      CompressionCodec codec = getCodec();
       OutputStream bos1 = null;
       if (downStreamBufferSize > 0) {
         bos1 = new BufferedOutputStream(downStream, downStreamBufferSize);
@@ -184,7 +182,8 @@
public final class Compression {
       else {
         bos1 = downStream;
       }
-      codec.getConf().setInt("io.file.buffer.size", 32 * 1024);
+      Configurable c = (Configurable) codec;
+      c.getConf().setInt("io.file.buffer.size", 32 * 1024);
       CompressionOutputStream cos =
           codec.createOutputStream(bos1, compressor);
       BufferedOutputStream bos2 =
